7da8f3
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -122,21 +122,25 @@
public static boolean isDedicatedCluster(Configuration conf) {
 
   public static SparkSession getSparkSession(HiveConf conf,
       SparkSessionManager sparkSessionManager) throws HiveException {
-    SparkSession sparkSession = SessionState.get().getSparkSession();
-    HiveConf sessionConf = SessionState.get().getConf();
 
-    // Spark configurations are updated close the existing session
-    // In case of async queries or confOverlay is not empty,
-    // sessionConf and conf are different objects
-    if (sessionConf.getSparkConfigUpdated() || conf.getSparkConfigUpdated()) {
-      sparkSessionManager.closeSession(sparkSession);
-      sparkSession = null;
-      conf.setSparkConfigUpdated(false);
-      sessionConf.setSparkConfigUpdated(false);
+    SessionState sessionState = SessionState.get();
+    synchronized (sessionState) {
+      SparkSession sparkSession = sessionState.getSparkSession();
+      HiveConf sessionConf = sessionState.getConf();
+
+      // Spark configurations are updated close the existing session
+      // In case of async queries or confOverlay is not empty,
+      // sessionConf and conf are different objects
+      if (sessionConf.getSparkConfigUpdated() || conf.getSparkConfigUpdated()) {
+        sparkSessionManager.closeSession(sparkSession);
+        sparkSession = null;
+        conf.setSparkConfigUpdated(false);
+        sessionConf.setSparkConfigUpdated(false);
+      }
+      sparkSession = sparkSessionManager.getSession(sparkSession, conf, true);
+      sessionState.setSparkSession(sparkSession);
+      return sparkSession;
     }
-    sparkSession = sparkSessionManager.getSession(sparkSession, conf, true);
-    SessionState.get().setSparkSession(sparkSession);
-    return sparkSession;
   }
 
   /**
